[fix][broker] Correct two race conditions in the tracker code and logic bug in InMemoryDelayedDeliveryTracker that failed with NoSuchElementException#25681
Conversation
39814a6 to
c11c32f
Compare
992ab28 to
0038cf4
Compare
…rMultipleConsumers - apache#25617 DelayedDeliveryTracker is not thread safe, and any access to it must be done holding the object lock. There were five cases I found, four of them I could correct just by adding synchronized to the method declaration. In one of them I used a manual scope since it was only a subset of the method. This is a significant issue, as exceptions here were uncaught and put the broker into an invalid state that prevented some delayed messages from being delivered until restart. The included unit test failed 100% of the time when run locally without the fix.
…Tracker While reviewing the code for more issues to explain the crashes we were still seeing, I noticed that InMemoryDelayedDeliveryTracker contains two major state fields: - delayedMessageMap - delayedMessagesCount These are kept in sync in various methods, but those methods were _not_ correctly synchronized, which meant it was trivial for us to get them out of sync and then crash peeking at the map. Added a unit test that triggers 100% of the time before the fix.
…o InMemoryDelayedDeliveryTracker
When the same ledger/entry messages come into InMemoryDelayedDeliveryTracker, we can desync the
delayedMessagesCount/delayedMessageMap count such that there are less items in the map than the count
If then we pop down to zero (not clear), we can get:
```
java.util.NoSuchElementException
at java.base/java.util.TreeMap.key(TreeMap.java:1637)
at java.base/java.util.TreeMap.firstKey(TreeMap.java:302)
at org.apache.pulsar.broker.delayed.InMemoryDelayedDeliveryTracker.nextDeliveryTime(InMemoryDelayedDeliveryTracker.java:291)
at org.apache.pulsar.broker.delayed.AbstractDelayedDeliveryTracker.updateTimer(AbstractDelayedDeliveryTracker.java:117)
at org.apache.pulsar.broker.delayed.InMemoryDelayedDeliveryTracker.getScheduledMessages(InMemoryDelayedDeliveryTracker.java:247)
at org.apache.pulsar.broker.delayed.InMemoryDeliveryTrackerTest.testAddMultipleMessagesSameWindow(InMemoryDeliveryTrackerTest.java:328)
```
Instead, we track if the bitmap doesn't already contain the item, and only increment delayedMessagesCount if not.
0038cf4 to
66df8a9
Compare
…tipleConsumersClassic with test
…DeliveryTracker" This reverts commit 522604d.
|
I have resolved all current PR review comments and am ready for a re-review. |
|
Thanks @lhotari - Do we have any idea when this fix might make it into a public release? Could it make it into a 4.x branch? We'd like to make some plans, and knowing what timetable to expect would be really helpful. |
|
Yes, it's a bug fix and it will be backported into 4.0.x (prob 4.0.11) release |
|
@chamons I have cherry-picked in this to branch-4.0, branch-4.1 and branch-4.2 . Since the implementations in each branch is slightly different, the change has been backported. Please note that 4.1.x is no longer supported, so the release won't be made on that branch although the fix is present on the branch. It's recommended to upgrade to 4.2.2 once it comes out within about a month. In the mean time, you can build your own release. The Pulsar release is performed using these steps: https://pulsar.apache.org/contribute/release-process/#build-release-artifacts . The "Build release artifacts" and "Stage docker images" are the only required steps in this case. |
Fixes #25617
Motivation
While running a production system under load, we found a traffic pattern that could cause uncaught exceptions in the brokers of our pulsar cluster. If you have a large spike of messages with a matching deliver_at_time, it is possible to trigger java.util.NoSuchElementException and other exceptions:
This is a significant issue, as exceptions here were uncaught and put the broker into an invalid state that prevented some delayed messages from being delivered until restart.
Using our external repro as a guide, we were able to isolate two race conditions and a logic bug that triggered exceptions that matched these.
With these three fixes, we were able to run our sample case for 30 minutes straight without exceptions (before hand it would trigger within 3-5 minutes).
Modifications
Posting each commit's description here for simplicity:
Verifying this change
I have a clean CI here
The included unit tests fail 100% of the time when run locally without the fix, and we were able to run https://github.com/chamons/pulsar-scheduled-exception-repro without issues with the fixes included.
Does this pull request potentially affect one of the following parts:
If the box was checked, please highlight the changes